[fix] Write stuck due to pending add callback by multiple threads#4557
[fix] Write stuck due to pending add callback by multiple threads#4557StevenLuMT merged 3 commits intoapache:masterfrom
Conversation
|
Great analysis @poorbarcode. IIRC, there might be some other pending PRs in this area where the review isn't complete. I'll check if they are related. |
|
There are a few open Issues/PRs that haven't been processed and which could be related:
@poorbarcode would you be able to check if they are related and relevant? |
|
rerun failure checks |
|
@poorbarcode please take a look at the tests - there failures/timeouts |
|
reopen's reason: rerun failure checks |
https://github.com/apache/bookkeeper/actions/runs/14370207115/job/40291989272?pr=4557 |
6be195f to
1632cab
Compare
|
Rebased master |
…ache#4557) [fix] Write stuck due to pending add callback by multiple threads (apache#4557) (cherry picked from commit e47926b) (cherry picked from commit 143e5a8)
…ache#4557) [fix] Write stuck due to pending add callback by multiple threads (apache#4557) (cherry picked from commit e47926b) (cherry picked from commit 143e5a8)

Motivation
Background: the normal steps of adding an entry
PendingAddOp.writeCompleteafter receiving the response from BK servers.Background: the steps of disconnection
PendingAddOp.writeComplete. You can reproduce this flow by the new testtestAddEntriesCallbackWithBKClientThreadIssue-1: write stuck due to pending add callback by multiple threads
332client->BK1client->BK2client-> BK3ack:1/3ack:2/3completesince ack quorum is2/3PendingAddOp.writeCompletethread:bookkeeper workersthread:client-server ioSince there are multiple threads that will trigger all successful callbacks in the pending queue, it may cause the following race condition[code-2]
thread-1andthread-2may be triggered by differentPendingAddOpsthread-1thread-2successsuccessqueue.popqueue.pop[1] code link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L307
[2] code-link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2092-L2124
Issue-2: ledger will be closed with a incorrect length
Since the task that triggers all successful callbacks in the pending queue may be run in
IOthread, the task "triggers all successful callbacks in the pending queue" and closing ledger may concurrectly executeworker-threadio-threadsuccessledger.lengthwhich was popped out from the queue[code-3]queue.popand pop nothingledger.LACThe variables
ledger.LACandledger.lengthdo not match[3] code-link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2076-L2084
The issue we encountered
A pulsar topic is stuck at
ClosingLedgerstatepulsar topic stats
{ "entriesAddedCounter" : 12485917, "numberOfEntries" : 126383, "totalSize" : 53668291, "currentLedgerEntries" : 137418, "currentLedgerSize" : 58374388, "lastLedgerCreatedTimestamp" : "2025-02-06T09:13:45.371Z", "waitingCursorsCount" : 1, "pendingAddEntriesCount" : 8342123, "lastConfirmedEntry" : "41901:126416", "state" : "ClosingLedger", "ledgers" : [ { "ledgerId" : 41901, "entries" : 0, "size" : 0, "offloaded" : false, "underReplicated" : false } ], "cursors" : { ...logs
Changes
Switch the thread to
Bookkeeper worksif the connection is broken.